f9d1b6
@@ -20,10 +20,14 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -79,7 +83,7 @@
public Path open(String user, Configuration conf)
       this.user = user;
       File t = File.createTempFile("templeton", null);
       tokenPath = new Path(t.toURI());
-      Token fsToken = getFSDelegationToken(user, conf);
+      Token[] fsToken = getFSDelegationToken(user, conf);
       String hcatTokenStr;
       try {
         hcatTokenStr = buildHcatDelegationToken(user);
@@ -130,11 +134,11 @@
public void addArgs(List<String> args) {
     }
   }
 
-  class TokenWrapper {
-    Token<?> token;
+  private static class TokenWrapper {
+    Token<?>[] tokens = new Token<?>[0];
   }
 
-  private Token<?> getFSDelegationToken(String user,
+  private Token<?>[] getFSDelegationToken(String user,
                       final Configuration conf)
     throws IOException, InterruptedException {
     LOG.info("user: " + user + " loginUser: " + UserGroupInformation.getLoginUser().getUserName());
@@ -142,18 +146,32 @@
public void addArgs(List<String> args) {
 
     final TokenWrapper twrapper = new TokenWrapper();
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
-      public Object run() throws IOException {
-        FileSystem fs = FileSystem.get(conf);
-        //todo: according to JavaDoc this seems like private API: addDelegationToken should be used
-        twrapper.token = fs.getDelegationToken(ugi.getShortUserName());
+      public Object run() throws IOException, URISyntaxException {
+        Credentials creds = new Credentials();
+        //get Tokens for default FS.  Not all FSs support delegation tokens, e.g. WASB
+        collectTokens(FileSystem.get(conf), twrapper, creds, ugi.getShortUserName());
+        //get tokens for all other known FSs since Hive tables may result in different ones
+        //passing "creds" prevents duplicate tokens from being added
+        Collection<String> URIs = conf.getStringCollection("mapreduce.job.hdfs-servers");
+        for(String uri : URIs) {
+          LOG.debug("Getting tokens for " + uri);
+          collectTokens(FileSystem.get(new URI(uri), conf), twrapper, creds, ugi.getShortUserName());
+        }
         return null;
       }
     });
-    return twrapper.token;
-
+    return twrapper.tokens;
   }
-
-  private void writeProxyDelegationTokens(final Token<?> fsToken,
+  private static void collectTokens(FileSystem fs, TokenWrapper twrapper, Credentials creds, String userName) throws IOException {
+    Token[] tokens = fs.addDelegationTokens(userName, creds);
+    if(tokens != null && tokens.length > 0) {
+      twrapper.tokens = ArrayUtils.addAll(twrapper.tokens, tokens);
+    }
+  }
+  /**
+   * @param fsTokens not null
+   */
+  private void writeProxyDelegationTokens(final Token<?> fsTokens[],
                       final Token<?> msToken,
                       final Configuration conf,
                       String user,
@@ -168,7 +186,9 @@
private void writeProxyDelegationTokens(final Token<?> fsToken,
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
       public Object run() throws IOException {
         Credentials cred = new Credentials();
-        cred.addToken(fsToken.getService(), fsToken);
+        for(Token<?> fsToken : fsTokens) {
+          cred.addToken(fsToken.getService(), fsToken);
+        }
         cred.addToken(msToken.getService(), msToken);
         cred.writeTokenStorageFile(tokenPath, conf);
         return null;
